d6f230
@@ -42,6 +42,7 @@
import org.apache.hadoop.hbase.zookeeper.ZKUtil;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
 import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.AuthFailedException;
 import org.apache.zookeeper.KeeperException.ConnectionLossException;
 import org.apache.zookeeper.KeeperException.SessionExpiredException;
 
@@ -226,6 +227,9 @@
public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     try {
       addresses = fetchSlavesAddresses(peer.getZkw());
     } catch (KeeperException ke) {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Fetch salves addresses failed.", ke);
+      }
       reconnectPeer(ke, peer);
       addresses = Collections.emptyList();
     }
@@ -332,7 +336,8 @@
public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
    * @param peer
    */
   private void reconnectPeer(KeeperException ke, ReplicationPeer peer) {
-    if (ke instanceof ConnectionLossException || ke instanceof SessionExpiredException) {
+    if (ke instanceof ConnectionLossException || ke instanceof SessionExpiredException
+        || ke instanceof AuthFailedException) {
       LOG.warn("Lost the ZooKeeper connection for peer " + peer.getClusterKey(), ke);
       try {
         peer.reloadZkWatcher();
@@ -491,4 +496,4 @@
public class ReplicationPeersZKImpl extends ReplicationStateZKBase implements Re
     }
 
   }
-}
\ No newline at end of file
+}
